Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Enable configuration of a CDC mutation info Callable for CDC Writes into BigQuery #32878

Open
wants to merge 15 commits into
base: master
Choose a base branch
from

Conversation

prodriguezdefino
Copy link
Contributor

@prodriguezdefino prodriguezdefino commented Oct 18, 2024

This change enables the configuration of a Callable function to populate the CDC mutation information on a per record basis when using BigQuery as destination with CDC writes.

Given that beam.io.WriteToBigQuery supports input data as Dict or beam.Row, the configured CDC info Callable should conform to one of those 2 types: Callable[[Dict], Dict] or Callable[[beam.Row], beam.Row].

Example configuration for Dict type data:

    ...
    data = [
        # record: (name, value)
        {
            'name': 'cdc_test', 'value': 5
        },
        {
            'name': 'cdc_test', 'value': 3
        }
    ]

    schema = {
        "fields": [{
            "name": "name", "type": "STRING"
        }, {
            "name": "value", "type": "INTEGER"
        }]
    }

    def cdc_info_fn(data: Dict) -> Dict:
      return {
          'mutation_type': 'UPSERT',
          'change_sequence_number': 'AAA/' + str(data["value"])
      }
    
    with beam.Pipeline(argv=self.args) as p:
      _ = (
          p
          | beam.Create(data)
          | beam.io.WriteToBigQuery(
              table=table_id,
              method=beam.io.WriteToBigQuery.Method.STORAGE_WRITE_API,
              use_cdc_writes=cdc_info_fn,
              schema=schema,
              primary_key=["name"],
              use_at_least_once=True))

# expected data written on BQ table after pipeline finishes
# {
#    'name': 'cdc_test', 'value': 5
# }

Analogously, in case of processing Beam Rows the pipeline would look like:

    ...
    row_data = [
        beam.Row(name="cdc_test", value=5, route=3),
        beam.Row(name="cdc_test", value=3, route=3),
        beam.Row(name="cdc_test", value=2, route=1)
    ]
    def cdc_info_rows(row: beam.Row) -> beam.Row:
      return beam.Row(
          mutation_type="UPSERT",
          change_sequence_number="AAA/" + str(row.value + row.route))

    with beam.Pipeline(argv=self.args) as p:
       _ = (
          p
          | beam.Create(row_data)
          | beam.io.WriteToBigQuery(
              table=table_id,
              method=beam.io.WriteToBigQuery.Method.STORAGE_WRITE_API,
              use_cdc_writes=cdc_info_rows,
              primary_key=["name"],
              use_at_least_once=True))

# expected data written on BQ table after pipeline finishes
# {
#    'name': 'cdc_test', 'value': 5, 'route': 3
# }

GitHub Actions Tests Status (on master branch)

Build python source distribution and wheels
Python tests
Java tests
Go tests

See CI.md for more information about GitHub Actions CI or the workflows README to see a list of phrases to trigger workflows.

@github-actions github-actions bot added the build label Oct 21, 2024
@prodriguezdefino prodriguezdefino changed the title [WIP] Enable configuration of mutation info callable for CDC Writes into BigQuery Enable configuration of mutation info callable for CDC Writes into BigQuery Oct 22, 2024
@prodriguezdefino prodriguezdefino changed the title Enable configuration of mutation info callable for CDC Writes into BigQuery Enable configuration of a CDC mutation info Callable for CDC Writes into BigQuery Oct 22, 2024
@prodriguezdefino prodriguezdefino marked this pull request as ready for review November 12, 2024 22:31
Copy link
Contributor

Assigning reviewers. If you would like to opt out of this review, comment assign to next reviewer:

R: @shunping for label python.
R: @Abacn for label build.
R: @johnjcasey for label io.

Available commands:

  • stop reviewer notifications - opt out of the automated review tooling
  • remind me after tests pass - tag the comment author after tests pass
  • waiting on author - shift the attention set back to the author (any comment or push by the author will return the attention set to the reviewers)

The PR bot will only process comments in the main thread (not review comments).

Copy link
Contributor

Reminder, please take a look at this pr: @shunping @Abacn @johnjcasey

Copy link
Contributor

Assigning new set of reviewers because Pr has gone too long without review. If you would like to opt out of this review, comment assign to next reviewer:

R: @jrmccluskey for label python.
R: @damccorm for label build.
R: @ahmedabu98 for label io.

Available commands:

  • stop reviewer notifications - opt out of the automated review tooling
  • remind me after tests pass - tag the comment author after tests pass
  • waiting on author - shift the attention set back to the author (any comment or push by the author will return the attention set to the reviewers)

Copy link
Contributor

github-actions bot commented Dec 2, 2024

Reminder, please take a look at this pr: @jrmccluskey @damccorm @ahmedabu98

@@ -2550,7 +2577,7 @@ def __init__(
use_at_least_once=False,
with_auto_sharding=False,
num_storage_api_streams=0,
use_cdc_writes: bool = False,
use_cdc_writes: UseCdcWrites = False,
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Could you add some more context on why we need this PR? What use case is it enabling which previously was impossible/is there a reason having this as a parameter is better than just recommending users do preprocessing before the write step? As much as we can, I'd prefer to keep custom user code out of the IO for debugability.

I'll also note that this parameter is now very overloaded. I'd probably prefer an extra param here over repurposing this one.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

But lets start with the first question, because I'm not sure about the change as a whole yet

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Originally this change was part of #32529 and given the push to get it included for the v2.60.0 release @ahmedabu98 recommended partition it in two different PRs.

Before #32529 Python SDK was not capable of using CDC writes into BQ and this change brings parity with BigQueryIO by exposing a lambda to include Row Mutation information as part of the row to be ingested. Currently, a SDK user would need to know how to structure their Rows to be ingested using CDC or, if they use Dicts as their data format, their provided schema should include the row mutation information making it not matching with the actual BigQuery table schema they want to write to (otherwise the xlang protocol wouldn't work).

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I'll also note that this parameter is now very overloaded. I'd probably prefer an extra param here over repurposing this one.

We have similar overloads in this same file, see here and also we have other instances on where the overload is silently added on a typeless argument.

The strict type definition simplifies the understanding of the shape for the argument, and to some degree the validations made later (would have been even simpler and more strict with Python > 3.10).

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Currently, a SDK user would need to know how to structure their Rows to be ingested using CDC or, if they use Dicts as their data format, their provided schema should include the row mutation information making it not matching with the actual BigQuery table schema they want to write to (otherwise the xlang protocol wouldn't work).

Isn't this still true with your change? The only difference is that they're now encapsulating the conversion logic in their Write transform instead of having a step to do it beforehand. Basically, as I understand it, the following 2 blocks are equivalent (and not particularly different to write):

pcoll
| WriteToBigQuery(..., use_cdc_writes=my_fn, ...)

and:

pcoll
| beam.Map(my_fn)
| WriteToBigQuery(..., use_cdc_writes=True, ...)

the main difference is that the 2nd is easier to debug IMO. Am I right that those are equivalent in all cases or am I missing something?

this change brings parity with BigQueryIO

This is a good reason to consider the change, but not enough to add it IMO

We have similar overloads in this same file, see here and also we have other instances on where the overload is silently added on a typeless argument.

To be honest, I don't love that usage either. However, there are a few differences there:

  1. It is not a user facing API
  2. table is a more general arg than use_cdc_writes (which is no longer descriptive with this change)

Copy link
Contributor

github-actions bot commented Dec 5, 2024

Assigning new set of reviewers because Pr has gone too long without review. If you would like to opt out of this review, comment assign to next reviewer:

R: @liferoad for label python.
R: @Abacn for label build.
R: @chamikaramj for label io.

Available commands:

  • stop reviewer notifications - opt out of the automated review tooling
  • remind me after tests pass - tag the comment author after tests pass
  • waiting on author - shift the attention set back to the author (any comment or push by the author will return the attention set to the reviewers)

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Projects
None yet
Development

Successfully merging this pull request may close these issues.

2 participants